Skip to content

Conversation

@Nikita-Shupletsov
Copy link
Contributor

@Nikita-Shupletsov Nikita-Shupletsov commented Jan 27, 2026

This PR fixes a bug when KS doesn't close stores if the shutdown was
triggered during rebalance where an active tasks gets converted to a
standby one and put into pendingTasksToInit

  • Added logic to close pending tasks to init.
  • Made standby task closure similar to the one for active tasks.
  • Added a separate method for getting standby tasks from task registry.
  • Added an integration test that reproduces the issue.

Reviewers: Matthias J. Sax matthias@confluent.io

Added logic to close pending tasks to init.
Made standby task closure similar to the one for active tasks.
Added a separate method for getting standby tasks from task registry.
Added an integration test that reproduces the issue.
@github-actions github-actions bot added triage PRs from the community streams labels Jan 27, 2026
@mjsax mjsax added ci-approved and removed triage PRs from the community labels Jan 29, 2026
Copy link
Member

@mjsax mjsax left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Made a first pass.

Can you update the PR description adding context on what the bug exactly is, and when we hit it? It seems to be related to not closing "pending tasks", but might be good to give some more context.

Nikita-Shupletsov and others added 2 commits January 29, 2026 21:52
…ms/integration/RebalanceTaskClosureIntegrationTest.java

Co-authored-by: Matthias J. Sax <mjsax@apache.org>
Copy link
Member

@mjsax mjsax left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Some more follow ups -- also need to get input from @lucasbru...

} else {
standbyTasks.add(pendingTask);
}
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for updating the PR description. It say "shutdown during rebalance when active task become standby tasks" but seems it goes either way, and the PR is actually fixing both direction?

Copy link
Member

@lucasbru lucasbru left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good find, thanks for identifying the problem.

An alternative way to fix it would be, I think, to consider pendingTasksToInit as a subset of Tasks.allTasks. Then conceptually, it may be simpler to say "close all tasks". But we'd have to check carefully what other places Tasks.allTasks and related methods are used, and if we are changing one of those places. In that case, we'd still need the "remove" fix you did.

@lucasbru
Copy link
Member

lucasbru commented Feb 3, 2026

An alternative way to fix it would be, I think, to consider pendingTasksToInit as a subset of Tasks.allTasks. Then conceptually, it may be simpler to say "close all tasks". But we'd have to check carefully what other places Tasks.allTasks and related methods are used, and if we are changing one of those places. In that case, we'd still need the "remove" fix you did.

On second thought, that would probably create more problems than it solves. But I wonder if we should rename "allTasks" to "allInitializedTasks" then?

@Nikita-Shupletsov
Copy link
Contributor Author

Using drainPendingTasksToInit sounds like a good idea to me. Not sure if there are any complications though, but it would seem cleaner code wise.

the problem with drain is that we always try to remove the task we are closing from the task registry: https://github.com/Nikita-Shupletsov/kafka/blob/8ee99ee82df2cf89fbb769d26c66395fd3a63761/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java#L1405

if we drain the pending tasks, there will be no way for us to confirm it ever existed there. so we either need to relax the check in the remove method or have a separate branch for closing pending to init tasks

@mjsax
Copy link
Member

mjsax commented Feb 3, 2026

maybe we would need to add a new member to TasksRegistry taking "pending tasks" which got closed?

Yes, that is what I mean by this -- after we drain the pending-tasks-to-init, we would add them into some new "pending-task-to-close" member inside TaskRegistry and update removeTask so check ownership based on this new member variable.

It's to some extend "cosmetics" as we only move all "pending task" from one collection into a different one, but it preserves the invariant we have, and makes the code cleaner.

@Nikita-Shupletsov
Copy link
Contributor Author

But I wonder if we should rename "allTasks" to "allInitializedTasks" then?
@lucasbru would you mind if I did that in a followup PR?

@lucasbru
Copy link
Member

lucasbru commented Feb 4, 2026

But I wonder if we should rename "allTasks" to "allInitializedTasks" then?
@lucasbru would you mind if I did that in a followup PR?

No problem, it's a minor thing.

Copy link
Member

@mjsax mjsax left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Few follow up comments. -- Still working through the integration test, but might be good to share what I have right away.

Nikita-Shupletsov and others added 4 commits February 5, 2026 14:48
Co-authored-by: Matthias J. Sax <mjsax@apache.org>
Co-authored-by: Matthias J. Sax <mjsax@apache.org>
Co-authored-by: Matthias J. Sax <mjsax@apache.org>
Modified the test so it actually reproduces the issue.
Small refactoring.
Nikita-Shupletsov and others added 6 commits February 5, 2026 20:37
Co-authored-by: Matthias J. Sax <mjsax@apache.org>
Co-authored-by: Matthias J. Sax <mjsax@apache.org>
…ms/integration/RebalanceTaskClosureIntegrationTest.java

Co-authored-by: Matthias J. Sax <mjsax@apache.org>
…ms/integration/RebalanceTaskClosureIntegrationTest.java

Co-authored-by: Matthias J. Sax <mjsax@apache.org>
…ms/integration/RebalanceTaskClosureIntegrationTest.java

Co-authored-by: Matthias J. Sax <mjsax@apache.org>
…ms/integration/RebalanceTaskClosureIntegrationTest.java

Co-authored-by: Matthias J. Sax <mjsax@apache.org>
…ms/integration/RebalanceTaskClosureIntegrationTest.java

Co-authored-by: Matthias J. Sax <mjsax@apache.org>
@mjsax mjsax merged commit 127a644 into apache:trunk Feb 6, 2026
23 checks passed
@mjsax
Copy link
Member

mjsax commented Feb 6, 2026

Thanks for the fix. -- Merged to trunk.

Cherry-picking to older branches is not clean. Can you maybe look into it and open new PRs all the way back to 4.2, 4.1, and 4.0 branches?

Nikita-Shupletsov added a commit to Nikita-Shupletsov/kafka that referenced this pull request Feb 6, 2026
This PR fixes a bug when KS doesn't close stores if the shutdown was
triggered during rebalance where an active tasks gets converted to a
standby one and put into pendingTasksToInit

* Added logic to close pending tasks to init.
* Made standby task closure similar to the one for active tasks.
* Added a separate method for getting standby tasks from task registry.
* Added an integration test that reproduces the issue.

Reviewers: Matthias J. Sax <matthias@confluent.io>

---------

Co-authored-by: Matthias J. Sax <mjsax@apache.org>
Nikita-Shupletsov added a commit to Nikita-Shupletsov/kafka that referenced this pull request Feb 6, 2026
This PR fixes a bug when KS doesn't close stores if the shutdown was
triggered during rebalance where an active tasks gets converted to a
standby one and put into pendingTasksToInit

* Added logic to close pending tasks to init.
* Made standby task closure similar to the one for active tasks.
* Added a separate method for getting standby tasks from task registry.
* Added an integration test that reproduces the issue.

Reviewers: Matthias J. Sax <matthias@confluent.io>

---------

Co-authored-by: Matthias J. Sax <mjsax@apache.org>
 Conflicts:
	streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java
Nikita-Shupletsov added a commit to Nikita-Shupletsov/kafka that referenced this pull request Feb 7, 2026
This PR fixes a bug when KS doesn't close stores if the shutdown was
triggered during rebalance where an active tasks gets converted to a
standby one and put into pendingTasksToInit

* Added logic to close pending tasks to init.
* Made standby task closure similar to the one for active tasks.
* Added a separate method for getting standby tasks from task registry.
* Added an integration test that reproduces the issue.

Reviewers: Matthias J. Sax <matthias@confluent.io>

---------

Co-authored-by: Matthias J. Sax <mjsax@apache.org>
 Conflicts:
	streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java
Nikita-Shupletsov added a commit to Nikita-Shupletsov/kafka that referenced this pull request Feb 7, 2026
This PR fixes a bug when KS doesn't close stores if the shutdown was
triggered during rebalance where an active tasks gets converted to a
standby one and put into pendingTasksToInit

* Added logic to close pending tasks to init.
* Made standby task closure similar to the one for active tasks.
* Added a separate method for getting standby tasks from task registry.
* Added an integration test that reproduces the issue.

Reviewers: Matthias J. Sax <matthias@confluent.io>

---------

Co-authored-by: Matthias J. Sax <mjsax@apache.org>
 Conflicts:
	streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java
@Nikita-Shupletsov
Copy link
Contributor Author

4.0: #21421
4.1: #21422
4.2: #21423

Nikita-Shupletsov added a commit to Nikita-Shupletsov/kafka that referenced this pull request Feb 9, 2026
This PR fixes a bug when KS doesn't close stores if the shutdown was
triggered during rebalance where an active tasks gets converted to a
standby one and put into pendingTasksToInit

* Added logic to close pending tasks to init.
* Made standby task closure similar to the one for active tasks.
* Added a separate method for getting standby tasks from task registry.
* Added an integration test that reproduces the issue.

Reviewers: Matthias J. Sax <matthias@confluent.io>

---------

Co-authored-by: Matthias J. Sax <mjsax@apache.org>
 Conflicts:
	streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java
clolov pushed a commit that referenced this pull request Feb 9, 2026
This PR fixes a bug when KS doesn't close stores if the shutdown was
triggered during rebalance where an active tasks gets converted to a
standby one and put into pendingTasksToInit

* Added logic to close pending tasks to init.
* Made standby task closure similar to the one for active tasks.
* Added a separate method for getting standby tasks from task registry.
* Added an integration test that reproduces the issue.

Reviewers: Matthias J. Sax <matthias@confluent.io>
Conflicts:

streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java
@Nikita-Shupletsov
Copy link
Contributor Author

@lucasbru, @mjsax the follow-up PR to rename the methods in TaskRegistry: #21448
thanks in advance!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants